76a3d5
@@ -2680,19 +2680,24 @@
private static void copyFiles(final HiveConf conf, final FileSystem destFs,
           name = itemName;
           filetype = "";
         }
-
-        final String srcGroup = srcFile.getGroup();
+        final boolean renameNonLocal = !needToCopy && !isSrcLocal;
+        // If we do a rename for a non-local file, we will be transfering the original
+        // file permissions from source to the destination. Else, in case of mvFile() where we
+        // copy from source to destination, we will inherit the destination's parent group ownership.
+        final String srcGroup = renameNonLocal ? srcFile.getGroup() :
+          fullDestStatus.getFileStatus().getGroup();
         if (null == pool) {
           Path destPath = new Path(destf, srcP.getName());
           try {
 
-            if (!needToCopy && !isSrcLocal) {
+            if (renameNonLocal) {
               for (int counter = 1; !destFs.rename(srcP,destPath); counter++) {
                 destPath = new Path(destf, name + ("_copy_" + counter) + filetype);
               }
             } else {
               destPath = mvFile(conf, srcP, destPath, isSrcLocal, srcFs, destFs, name, filetype);
             }
+
             if (null != newFiles) {
               newFiles.add(destPath);
             }
@@ -2706,7 +2711,7 @@
private static void copyFiles(final HiveConf conf, final FileSystem destFs,
             public ObjectPair<Path, Path> call() throws Exception {
               SessionState.setCurrentSessionState(parentSession);
               Path destPath = new Path(destf, srcP.getName());
-              if (!needToCopy && !isSrcLocal) {
+              if (renameNonLocal) {
                 for (int counter = 1; !destFs.rename(srcP,destPath); counter++) {
                   destPath = new Path(destf, name + ("_copy_" + counter) + filetype);
                 }
